d5a347
@@ -766,22 +766,27 @@
public class HConnectionManager {
         synchronized (this) {
           if (batchPool == null) {
             int maxThreads = conf.getInt("hbase.hconnection.threads.max", 256);
-            int coreThreads = conf.getInt("hbase.hconnection.threads.core", 0);
+            int coreThreads = conf.getInt("hbase.hconnection.threads.core", 256);
             if (maxThreads == 0) {
               maxThreads = Runtime.getRuntime().availableProcessors() * 8;
             }
-            long keepAliveTime = conf.getLong("hbase.hconnection.threads.keepalivetime", 10);
+            if (coreThreads == 0) {
+              coreThreads = Runtime.getRuntime().availableProcessors() * 8;
+            }
+            long keepAliveTime = conf.getLong("hbase.hconnection.threads.keepalivetime", 60);
             LinkedBlockingQueue<Runnable> workQueue =
               new LinkedBlockingQueue<Runnable>(maxThreads *
                 conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
                   HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS));
-            this.batchPool = new ThreadPoolExecutor(
+            ThreadPoolExecutor tpe = new ThreadPoolExecutor(
                 coreThreads,
                 maxThreads,
                 keepAliveTime,
                 TimeUnit.SECONDS,
                 workQueue,
                 Threads.newDaemonThreadFactory(toString() + "-shared-"));
+            tpe.allowCoreThreadTimeOut(true);
+            this.batchPool = tpe;
           }
           this.cleanupPool = true;
         }
